3672a2
@@ -32,7 +32,7 @@
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.hive.common.HiveStatsUtils;
 import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
@@ -46,7 +46,6 @@
 import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer.TableSpec;
 import org.apache.hadoop.hive.ql.plan.StatsNoJobWork;
 import org.apache.hadoop.hive.ql.plan.api.StageType;
-import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hadoop.mapred.FileSplit;
 import org.apache.hadoop.mapred.InputFormat;
 import org.apache.hadoop.mapred.InputSplit;
@@ -144,9 +143,8 @@
public void run() {
         long fileSize = 0;
         long numFiles = 0;
         FileSystem fs = dir.getFileSystem(conf);
-        List<FileStatus> fileList = 
-          ShimLoader.getHadoopShims().listLocatedStatus(fs, dir,
-                                                        hiddenFileFilter);
+        FileStatus[] fileList = HiveStatsUtils.getFileStatusRecurse(dir, -1, fs);
+
         boolean statsAvailable = false;
         for(FileStatus file: fileList) {
           if (!file.isDir()) {
@@ -155,7 +153,6 @@
public void run() {
             InputSplit dummySplit = new FileSplit(file.getPath(), 0, 0,
                 new String[] { partn.getLocation() });
             org.apache.hadoop.mapred.RecordReader<?, ?> recordReader =
-                (org.apache.hadoop.mapred.RecordReader<?, ?>)
                 inputFormat.getRecordReader(dummySplit, jc, Reporter.NULL);
             StatsProvidingRecordReader statsRR;
             if (recordReader instanceof StatsProvidingRecordReader) {
@@ -242,9 +239,8 @@
private int aggregateStats(ExecutorService threadPool) {
           long fileSize = 0;
           long numFiles = 0;
           FileSystem fs = dir.getFileSystem(conf);
-          List<FileStatus> fileList = 
-            ShimLoader.getHadoopShims().listLocatedStatus(fs, dir,
-                                                          hiddenFileFilter);
+          FileStatus[] fileList = HiveStatsUtils.getFileStatusRecurse(dir, -1, fs);
+
           boolean statsAvailable = false;
           for(FileStatus file: fileList) {
             if (!file.isDir()) {
@@ -252,8 +248,8 @@
private int aggregateStats(ExecutorService threadPool) {
                   table.getInputFormatClass(), jc);
               InputSplit dummySplit = new FileSplit(file.getPath(), 0, 0, new String[] { table
                   .getDataLocation().toString() });
-              org.apache.hadoop.mapred.RecordReader<?, ?> recordReader = (org.apache.hadoop.mapred.RecordReader<?, ?>) inputFormat
-                  .getRecordReader(dummySplit, jc, Reporter.NULL);
+              org.apache.hadoop.mapred.RecordReader<?, ?> recordReader =
+                  inputFormat.getRecordReader(dummySplit, jc, Reporter.NULL);
               StatsProvidingRecordReader statsRR;
               if (recordReader instanceof StatsProvidingRecordReader) {
                 statsRR = (StatsProvidingRecordReader) recordReader;
@@ -354,13 +350,6 @@
private void shutdownAndAwaitTermination(ExecutorService threadPool) {
     }
   }
 
-  private static final PathFilter hiddenFileFilter = new PathFilter() {
-    public boolean accept(Path p) {
-      String name = p.getName();
-      return !name.startsWith("_") && !name.startsWith(".");
-    }
-  };
-
   private String toString(Map<String, String> parameters) {
     StringBuilder builder = new StringBuilder();
     for (String statType : StatsSetupConst.supportedStats) {
